Skip to content

Conversation

@chenjian2664
Copy link
Contributor

@chenjian2664 chenjian2664 commented Apr 10, 2023

Description

#16709

Additional context and related issues

This pr is going to support MERGE for jdbc based connectors, follows the #16693. Here are some major changes made in this pr:
JdbcClient & BaseJdbcClient:

  1. Add getPrimaryKeys method to get primary keys for the merge target table, the method must not return empty if the connector declares supporting merge. The default implementation in BaseJdbcClient, and extracted function extractJdbcHandlesFromResultSet from getColumns for reuse it. Currently we use primary keys for perform delete and update for the merge.

  2. Using return result of getPrimaryKeys to check if the connector support merge.

  3. Add methods beginDeleteTableForMerge and finishDeleteTableForMerge for supporting FTE about the merge, the logic will be used if the connector supports retry and use transactional insert. The default implementation in BaseJdbcClient :

    1. beginDeleteTableForMerge method uses primary keys to build a temporary table

    2. finishDeleteTableForMerge uses syntax DELETE FROM merge_target WHERE EXISTS (SELECT 1 FROM (temp_table_data ) temp WHERE "getConjunctsBetweenTargetAndTemporaryTable(merge_target, temp)" ) to perform actual delete.

      temp_table_data is the sth like SELECT * FROM temp_table WHERE EXISTS (SELECT 1 FROM page_sink_table WHERE page_sink_table.id_column = temp_table.id_column) The logic is the same as original INSERT operation that using temporary table(INSERT INTO insert_target SELECT * FROM temp_table WHERE EXISTS (SELECT 1 FROM page_sink_table WHERE page_sink_table.id_column = temp_table.id_column)), the temp_table is built in beginDeleteTableForMerge

      The connectors returns special primary keys that can not build temporary table with get primary keys directly , i.e PostgreSql returns ctid as the primary key, it is a hidden column for every table, while building temporary table storing the column data we need to rename the special column for avoiding conflicts, in PostgreSql we build the temporary table with column name ctid_for_delete_merge to store the ctid values, then use getConjunctsBetweenTargetAndTemporaryTable to use correct column name to build the condition between target and temp table .

  4. Add method buildMergeRowIdConjuncts for the connectors that not support(or disable) building temporary table to perform delete, the method builds the WHERE condition for clause DELETE FROM merge_target WHERE... using primary keys (called in JdbcMergeSink )

  5. The method updateScanColumnsForMerge is the same as we support merge in Phoenix, it updates the scans for including the all the primary keys if possible.

DefaultJdbcMetadata:

  1. Add a deleteForMergeRollbackAction field for the rollback about the delete operation if the connectors supports delete using temporary table.
  2. The method getMergeRowIdColumnHandle will check if the connector support merge, if yes it will return the column handle that composed by the primary keys(returned by getPrimaryKeys in client)
  3. beginMerge will pass the all the infos that needed by the merge, it's similar to we do in Phoenix connector, the difference is we add a beginDeleteMerge process for connectors that will use temporary table to perform delete.
  4. finishMerge calls the finishInsert and finishDeleteForMerge

JdbcMergeSink:
It's moved from the Phoenix implementation, difference are:

  • the creating delete sink will return the insert sink with the temporary table handle if the connectors support delete using temporary table.
  • The update will use syntax UPDATE merge_target SET reserved_columns WHERE..., the reserved_columns are the columns who are not the primary keys, the values come from channels that provided by getReservedChannelsForUpdate , and the WHERE condition is built by jdbcClient.buildMergeRowIdConjuncts.

The connectors:
Implementation about the getPrimaryKeys :

  • PostgreSql using the ctid
  • Oracle using the ROWID
  • Ignite using the primary keys defined in table
  • Phoenix using the rowkey defined in table
    Other Jdbc connectors Like Mysql actual can perform the merge as long as the table has defined the primary key, but since it is not forced by the connector, we currently let the BaseJdbcClient.supportsMerge return false by default and not declare it supports merge.

Tests:
PostgreSql and Oracle support FTE, so the defined the supportMerge in the BaseJdbcFailureRecoveryTest for indicating the behavior, aslo update the test cases through the method in BaseJdbcFailureRecoveryTest.
Adjust lots of merge tests in Oracle due to naming length constraints.

Release notes

( ) This is not user-visible or docs only and no release notes are required.
(x) Release notes are required, please propose a release note for me.
( ) Release notes are required, with the following suggested text:

# Base Jdbc
* Support SQL MERGE for base jdbc connectors ({issue}`16709`)
* Support update/merge for PostgreSql
* Support update/merge for Ignite
* Support update/merge for Oracle
* Update Phoenix merge implementation by reusing base Jdbc merge implementation.

@cla-bot cla-bot bot added the cla-signed label Apr 10, 2023
@chenjian2664 chenjian2664 marked this pull request as draft April 10, 2023 10:52
@chenjian2664 chenjian2664 self-assigned this Apr 12, 2023
@github-actions github-actions bot added the docs label May 22, 2023
@chenjian2664 chenjian2664 marked this pull request as ready for review May 22, 2023 07:48
@kokosing
Copy link
Member

I was thinking to merge #16445 first. Is that ok to you?

@chenjian2664
Copy link
Contributor Author

I was thinking to merge #16445 first. Is that ok to you?

Sure!

@sajjoseph
Copy link
Contributor

@kokosing,
Now that #16445 is merged, shall we give this PR another try?

@chenjian2664
Obviously, SQL Server is missing from the list above.
Is there any reason why it was dropped other than it was just not implemented?

Thanks!

@kokosing
Copy link
Member

@chenjian2664 Would you like to rebase this PR?

I know that @vlad-lyutenko is going to improve UPDATE a bit still. There are some rough edges that we would like to improve.

@chenjian2664
Copy link
Contributor Author

@chenjian2664 Obviously, SQL Server is missing from the list above. Is there any reason why it was dropped other than it was just not implemented?

I remember it's because the performance is too bad on SQL Server

@chenjian2664
Copy link
Contributor Author

@chenjian2664 Would you like to rebase this PR?

Sure

I know that @vlad-lyutenko is going to improve UPDATE a bit still. There are some rough edges that we would like to improve.

Are you suggest that we could keep on suspend this PR?

@kokosing
Copy link
Member

Are you suggest that we could keep on suspend this PR?

It is was suspended already too long. It is independent effort.

@chenjian2664 chenjian2664 marked this pull request as draft October 18, 2023 15:58
@chenjian2664 chenjian2664 marked this pull request as ready for review November 17, 2023 09:39
Support update/merge for Postgresql
Support update/merge for Ignite
Support update/merge for Oracle
Refactor Phoenix merge implementation by reuse base jdbc merge implementation.
@github-actions
Copy link

This pull request has gone a while without any activity. Tagging the Trino developer relations team: @bitsondatadev @colebow @mosabua

@github-actions github-actions bot added the stale label Jan 16, 2024
@mosabua
Copy link
Member

mosabua commented Jan 16, 2024

👋 @kokosing @chenjian2664 @sajjoseph can you collaborate to figure out best next steps here?

@github-actions github-actions bot removed the stale label Jan 18, 2024
@chenjian2664 chenjian2664 mentioned this pull request Feb 1, 2024
@github-actions
Copy link

github-actions bot commented Feb 9, 2024

This pull request has gone a while without any activity. Tagging the Trino developer relations team: @bitsondatadev @colebow @mosabua

@github-actions github-actions bot added the stale label Feb 9, 2024
@github-actions
Copy link

github-actions bot commented Mar 1, 2024

Closing this pull request, as it has been stale for six weeks. Feel free to re-open at any time.

@github-actions github-actions bot closed this Mar 1, 2024
@wendigo wendigo reopened this Mar 1, 2024
@hashhar
Copy link
Member

hashhar commented Mar 4, 2024

@wendigo The work is continued at #20532

@wendigo wendigo closed this Mar 4, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Development

Successfully merging this pull request may close these issues.

6 participants